Skip to content

Conversation

wForget
Copy link
Member

@wForget wForget commented Sep 1, 2025

Which issue does this PR close?

Closes #2271.

Rationale for this change

Currently we prefer to use jvm-based libhdfs to implement native hdfs reader, which means we can support more hadoop file systems. But currently we hardcode to support only hdfs scheme, I want to make the supported hadoop file system schemes configurable.

What changes are included in this PR?

Make supported hadoop filesystem schemes configurable

How are these changes tested?

After patch #2244, the newly added test cases were successfully run

@codecov-commenter
Copy link

codecov-commenter commented Sep 1, 2025

Codecov Report

❌ Patch coverage is 37.50000% with 5 lines in your changes missing coverage. Please review.
✅ Project coverage is 57.90%. Comparing base (f09f8af) to head (d442a93).
⚠️ Report is 455 commits behind head on main.

Files with missing lines Patch % Lines
...la/org/apache/comet/objectstore/NativeConfig.scala 0.00% 5 Missing ⚠️
Additional details and impacted files
@@             Coverage Diff              @@
##               main    #2272      +/-   ##
============================================
+ Coverage     56.12%   57.90%   +1.77%     
- Complexity      976     1291     +315     
============================================
  Files           119      146      +27     
  Lines         11743    13376    +1633     
  Branches       2251     2374     +123     
============================================
+ Hits           6591     7745    +1154     
- Misses         4012     4374     +362     
- Partials       1140     1257     +117     

☔ View full report in Codecov by Sentry.
📢 Have feedback on the report? Share it here.

🚀 New features to boost your workflow:
  • ❄️ Test Analytics: Detect flaky tests, report on failures, and find test suite problems.

@wForget wForget marked this pull request as ready for review September 2, 2025 02:15
@comphead
Copy link
Contributor

comphead commented Sep 2, 2025

@parthchandra cc

}
.map_err(|e| ExecutionError::GeneralError(e.to_string()))?;
let (object_store, object_store_path): (Box<dyn ObjectStore>, Path) =
if is_hdfs_scheme(&url, object_store_configs) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is a little gotcha here when the scheme is s3a. In s3a's case, we replace s3a with s3 so that we can use the native object store implementation (

). If the user has s3a in the list of hdfs urls because they want to use the hadoop-aws implementation, then they will still end up with the native implementation.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, I made some adjustments, could you take another look?

@@ -40,6 +41,8 @@ object NativeConfig {
// Azure Data Lake Storage Gen2 secure configurations (can use both prefixes)
"abfss" -> Seq("fs.abfss.", "fs.abfs."))

val COMET_LIBHDFS_SCHEMES_KEY = "fs.comet.libhdfs.schemes"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we make this a Comet conf (i.e add it in CometConf so it is automatically documented)?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, moved to CometConf.

conf(s"spark.hadoop.$COMET_LIBHDFS_SCHEMES_KEY")
.doc(
"Defines filesystem schemes (e.g., hdfs, webhdfs) that the native side accesses " +
"via libhdfs, separated by commas.")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: perhaps we can mention that this configuration is valid only if comet has been built with the hdfs feature flag enabled.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, added this description

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Support additional hadoop file systems
4 participants